9eb25b
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.snapshot;
 
+import com.google.protobuf.CodedInputStream;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -30,7 +32,6 @@
import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -41,6 +42,7 @@
import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -68,7 +70,9 @@
import org.apache.hadoop.hbase.util.Threads;
 public final class SnapshotManifest {
   private static final Log LOG = LogFactory.getLog(SnapshotManifest.class);
 
-  private static final String DATA_MANIFEST_NAME = "data.manifest";
+  public static final String SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY = "snapshot.manifest.size.limit";
+
+  public static final String DATA_MANIFEST_NAME = "data.manifest";
 
   private List<SnapshotRegionManifest> regionManifests;
   private SnapshotDescription desc;
@@ -78,6 +82,7 @@
public final class SnapshotManifest {
   private final Configuration conf;
   private final Path workingDir;
   private final FileSystem fs;
+  private int manifestSizeLimit;
 
   private SnapshotManifest(final Configuration conf, final FileSystem fs,
       final Path workingDir, final SnapshotDescription desc,
@@ -87,6 +92,8 @@
public final class SnapshotManifest {
     this.workingDir = workingDir;
     this.conf = conf;
     this.fs = fs;
+
+    this.manifestSizeLimit = conf.getInt(SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 64 * 1024 * 1024);
   }
 
   /**
@@ -512,7 +519,9 @@
public final class SnapshotManifest {
     FSDataInputStream in = null;
     try {
       in = fs.open(new Path(workingDir, DATA_MANIFEST_NAME));
-      return SnapshotDataManifest.parseFrom(in);
+      CodedInputStream cin = CodedInputStream.newInstance(in);
+      cin.setSizeLimit(manifestSizeLimit);
+      return SnapshotDataManifest.parseFrom(cin);
     } catch (FileNotFoundException e) {
       return null;
     } finally {
